441b29
@@ -28,7 +28,6 @@
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -400,24 +399,31 @@
static String getPartitionName(Path tablePath, Path partitionPath,
    *          Specify how deep the search goes.
    * @throws IOException
    *           Thrown if we can't get lists from the fs.
-   * @throws HiveException 
+   * @throws HiveException
    */
 
   private void checkPartitionDirs(Path basePath, Set<Path> allDirs, int maxDepth) throws IOException, HiveException {
     ConcurrentLinkedQueue<Path> basePaths = new ConcurrentLinkedQueue<>();
     basePaths.add(basePath);
-    Set<Path> dirSet = Collections.newSetFromMap(new ConcurrentHashMap<Path, Boolean>());    
+    Set<Path> dirSet = Collections.newSetFromMap(new ConcurrentHashMap<Path, Boolean>());
     // Here we just reuse the THREAD_COUNT configuration for
     // HIVE_MOVE_FILES_THREAD_COUNT
-    final ExecutorService pool = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25) > 0 ? Executors
-        .newFixedThreadPool(conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25),
-            new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MSCK-GetPaths-%d").build())
-            : null;
+    int poolSize = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 15);
+
+    // Check if too low config is provided for move files. 2x CPU is reasonable max count.
+    poolSize = poolSize == 0 ? poolSize : Math.max(poolSize,
+        Runtime.getRuntime().availableProcessors() * 2);
+
+    // Fixed thread pool on need basis
+    final ThreadPoolExecutor pool = poolSize > 0 ? (ThreadPoolExecutor)
+        Executors.newFixedThreadPool(poolSize,
+            new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MSCK-GetPaths-%d").build()) : null;
+
     if (pool == null) {
       LOG.debug("Not-using threaded version of MSCK-GetPaths");
     } else {
       LOG.debug("Using threaded version of MSCK-GetPaths with number of threads "
-          + ((ThreadPoolExecutor) pool).getPoolSize());
+          + pool.getMaximumPoolSize());
     }
     checkPartitionDirs(pool, basePaths, dirSet, basePath.getFileSystem(conf), maxDepth, maxDepth);
     if (pool != null) {
@@ -427,11 +433,30 @@
private void checkPartitionDirs(Path basePath, Set<Path> allDirs, int maxDepth)
   }
 
   // process the basePaths in parallel and then the next level of basePaths
-  private void checkPartitionDirs(final ExecutorService pool,
+  private void checkPartitionDirs(final ThreadPoolExecutor pool,
       final ConcurrentLinkedQueue<Path> basePaths, final Set<Path> allDirs,
       final FileSystem fs, final int depth, final int maxDepth) throws IOException, HiveException {
     final ConcurrentLinkedQueue<Path> nextLevel = new ConcurrentLinkedQueue<>();
-    if (null == pool) {
+
+    // Check if thread pool can be used.
+    boolean useThreadPool = false;
+    if (pool != null) {
+      synchronized (pool) {
+        // In case of recursive calls, it is possible to deadlock with TP. Check TP usage here.
+        if (pool.getActiveCount() < pool.getMaximumPoolSize()) {
+          useThreadPool = true;
+        }
+
+        if (!useThreadPool) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Not using threadPool as active count:" + pool.getActiveCount()
+                + ", max:" + pool.getMaximumPoolSize());
+          }
+        }
+      }
+    }
+
+    if (null == pool || !useThreadPool) {
       for (final Path path : basePaths) {
         FileStatus[] statuses = fs.listStatus(path, FileUtils.HIDDEN_FILES_PATH_FILTER);
         boolean fileFound = false;
